Skip to content

Conversation

@kaviya2510
Copy link
Contributor

Added translation support for taskloop construct.

@kaviya2510 kaviya2510 marked this pull request as draft November 7, 2025 07:23
@llvmbot
Copy link
Member

llvmbot commented Nov 7, 2025

@llvm/pr-subscribers-flang-fir-hlfir
@llvm/pr-subscribers-mlir-openmp

@llvm/pr-subscribers-mlir-llvm

Author: Kaviya Rajendiran (kaviya2510)

Changes

Added translation support for taskloop construct.


Patch is 32.13 KiB, truncated to 20.00 KiB below, full version: https://github.com/llvm/llvm-project/pull/166903.diff

6 Files Affected:

  • (modified) llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h (+16)
  • (modified) llvm/include/llvm/Frontend/OpenMP/OMPKinds.def (+1)
  • (modified) llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp (+199)
  • (modified) mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp (+254-3)
  • (added) mlir/test/Target/LLVMIR/openmp-taskloop.mlir (+151)
  • (modified) mlir/test/Target/LLVMIR/openmp-todo.mlir (+1-14)
diff --git a/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h b/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
index b3d7ab4acf303..18828380abd32 100644
--- a/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
+++ b/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
@@ -1359,6 +1359,22 @@ class OpenMPIRBuilder {
         : DepKind(DepKind), DepValueType(DepValueType), DepVal(DepVal) {}
   };
 
+  /// Generator for `#omp taskloop`
+  ///
+  /// \param Loc The location where the taskloop construct was encountered.
+  /// \param AllocaIP The insertion point to be used for alloca instructions.
+  /// \param BodyGenCB Callback that will generate the region code.
+  /// \param LoopInfo Callback that return the CLI
+  /// \param LBVal Lowerbound value of loop
+  /// \param UBVal Upperbound value of loop
+  /// \param StepVal Step value of loop
+  /// \param Tied True if the task is tied, false if the task is untied.
+  LLVM_ABI InsertPointOrErrorTy createTaskloop(
+      const LocationDescription &Loc, InsertPointTy AllocaIP,
+      BodyGenCallbackTy BodyGenCB,
+      llvm::function_ref<llvm::Expected<llvm::CanonicalLoopInfo *>()> LoopInfo,
+      Value *LBVal, Value *UBVal, Value *StepVal, bool Tied = true);
+
   /// Generator for `#omp task`
   ///
   /// \param Loc The location where the task construct was encountered.
diff --git a/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def b/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
index 46b3d53a4b408..032495dfe9d61 100644
--- a/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
+++ b/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
@@ -95,6 +95,7 @@ __OMP_STRUCT_TYPE(KernelArgs, __tgt_kernel_arguments, false, Int32, Int32, VoidP
 __OMP_STRUCT_TYPE(AsyncInfo, __tgt_async_info, false, Int8Ptr)
 __OMP_STRUCT_TYPE(DependInfo, kmp_dep_info, false, SizeTy, SizeTy, Int8)
 __OMP_STRUCT_TYPE(Task, kmp_task_ompbuilder_t, false, VoidPtr, VoidPtr, Int32, VoidPtr, VoidPtr)
+__OMP_STRUCT_TYPE(Taskloop, kmp_task_info, false, VoidPtr, VoidPtr, Int32, VoidPtr, VoidPtr, Int64, Int64, Int64)
 __OMP_STRUCT_TYPE(ConfigurationEnvironment, ConfigurationEnvironmentTy, false,
                   Int8, Int8, Int8, Int32, Int32, Int32, Int32, Int32, Int32)
 __OMP_STRUCT_TYPE(DynamicEnvironment, DynamicEnvironmentTy, false, Int16)
diff --git a/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp b/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
index fff9a815e5368..e88e722b1370e 100644
--- a/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
+++ b/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
@@ -1933,6 +1933,205 @@ static Value *emitTaskDependencies(
   return DepArray;
 }
 
+OpenMPIRBuilder::InsertPointOrErrorTy OpenMPIRBuilder::createTaskloop(
+    const LocationDescription &Loc, InsertPointTy AllocaIP,
+    BodyGenCallbackTy BodyGenCB,
+    llvm::function_ref<llvm::Expected<llvm::CanonicalLoopInfo *>()> loopInfo,
+    Value *LBVal, Value *UBVal, Value *StepVal, bool Tied) {
+
+  if (!updateToLocation(Loc))
+    return InsertPointTy();
+
+  uint32_t SrcLocStrSize;
+  Constant *SrcLocStr = getOrCreateSrcLocStr(Loc, SrcLocStrSize);
+  Value *Ident = getOrCreateIdent(SrcLocStr, SrcLocStrSize);
+
+  BasicBlock *TaskloopExitBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.exit");
+  BasicBlock *TaskloopBodyBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.body");
+  BasicBlock *TaskloopAllocaBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.alloca");
+
+  InsertPointTy TaskloopAllocaIP =
+      InsertPointTy(TaskloopAllocaBB, TaskloopAllocaBB->begin());
+  InsertPointTy TaskloopBodyIP =
+      InsertPointTy(TaskloopBodyBB, TaskloopBodyBB->begin());
+
+  if (Error Err = BodyGenCB(TaskloopAllocaIP, TaskloopBodyIP))
+    return Err;
+
+  llvm::Expected<llvm::CanonicalLoopInfo *> result = loopInfo();
+  if (!result) {
+    return result.takeError();
+  }
+
+  llvm::CanonicalLoopInfo *CLI = result.get();
+  OutlineInfo OI;
+  OI.EntryBB = TaskloopAllocaBB;
+  OI.OuterAllocaBB = AllocaIP.getBlock();
+  OI.ExitBB = TaskloopExitBB;
+
+  // Add the thread ID argument.
+  SmallVector<Instruction *, 4> ToBeDeleted;
+  // dummy instruction to be used as a fake argument
+  OI.ExcludeArgsFromAggregate.push_back(createFakeIntVal(
+      Builder, AllocaIP, ToBeDeleted, TaskloopAllocaIP, "global.tid", false));
+
+  OI.PostOutlineCB = [this, Ident, LBVal, UBVal, StepVal, Tied,
+                      TaskloopAllocaBB, CLI, Loc,
+                      ToBeDeleted](Function &OutlinedFn) mutable {
+    // Replace the Stale CI by appropriate RTL function call.
+    assert(OutlinedFn.hasOneUse() &&
+           "there must be a single user for the outlined function");
+    CallInst *StaleCI = cast<CallInst>(OutlinedFn.user_back());
+
+    // HasShareds is true if any variables are captured in the outlined region,
+    // false otherwise.
+    bool HasShareds = StaleCI->arg_size() > 1;
+    Builder.SetInsertPoint(StaleCI);
+
+    // Gather the arguments for emitting the runtime call for
+    // @__kmpc_omp_task_alloc
+    Function *TaskAllocFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_omp_task_alloc);
+
+    Value *ThreadID = getOrCreateThreadID(Ident);
+
+    // Emit runtime call for @__kmpc_taskgroup
+    Function *TaskgroupFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_taskgroup);
+    Builder.CreateCall(TaskgroupFn, {Ident, ThreadID});
+
+    // The flags are set to 1 if the task is tied, 0 otherwise.
+    Value *Flags = Builder.getInt32(Tied);
+
+    Value *TaskSize = Builder.getInt64(
+        divideCeil(M.getDataLayout().getTypeSizeInBits(Taskloop), 8));
+
+    Value *SharedsSize = Builder.getInt64(0);
+    if (HasShareds) {
+      AllocaInst *ArgStructAlloca =
+          dyn_cast<AllocaInst>(StaleCI->getArgOperand(1));
+      assert(ArgStructAlloca &&
+             "Unable to find the alloca instruction corresponding to arguments "
+             "for extracted function");
+      StructType *ArgStructType =
+          dyn_cast<StructType>(ArgStructAlloca->getAllocatedType());
+      assert(ArgStructType && "Unable to find struct type corresponding to "
+                              "arguments for extracted function");
+      SharedsSize =
+          Builder.getInt64(M.getDataLayout().getTypeStoreSize(ArgStructType));
+    }
+
+    // Emit the @__kmpc_omp_task_alloc runtime call
+    // The runtime call returns a pointer to an area where the task captured
+    // variables must be copied before the task is run (TaskData)
+    CallInst *TaskData = Builder.CreateCall(
+        TaskAllocFn, {/*loc_ref=*/Ident, /*gtid=*/ThreadID, /*flags=*/Flags,
+                      /*sizeof_task=*/TaskSize, /*sizeof_shared=*/SharedsSize,
+                      /*task_func=*/&OutlinedFn});
+
+    // Get the pointer to loop lb, ub, step from task ptr
+    // and set up the lowerbound,upperbound and step values
+    llvm::Value *lb =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 5);
+    //  Value *LbVal_ext = Builder.CreateSExt(LBVal, Builder.getInt64Ty());
+    Builder.CreateStore(LBVal, lb);
+
+    llvm::Value *ub =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 6);
+    Builder.CreateStore(UBVal, ub);
+
+    llvm::Value *step =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 7);
+    Value *Step_ext = Builder.CreateSExt(StepVal, Builder.getInt64Ty());
+    Builder.CreateStore(Step_ext, step);
+    llvm::Value *loadstep = Builder.CreateLoad(Builder.getInt64Ty(), step);
+
+    if (HasShareds) {
+      Value *Shareds = StaleCI->getArgOperand(1);
+      Align Alignment = TaskData->getPointerAlignment(M.getDataLayout());
+      Value *TaskShareds = Builder.CreateLoad(VoidPtr, TaskData);
+      Builder.CreateMemCpy(TaskShareds, Alignment, Shareds, Alignment,
+                           SharedsSize);
+    }
+
+    // set up the arguments for emitting kmpc_taskloop runtime call
+    // setting default values for ifval, nogroup, sched, grainsize, task_dup
+    Value *IfVal = Builder.getInt32(1);
+    Value *NoGroup = Builder.getInt32(1);
+    Value *Sched = Builder.getInt32(0);
+    Value *GrainSize = Builder.getInt64(0);
+    Value *TaskDup = Constant::getNullValue(Builder.getPtrTy());
+
+    Value *Args[] = {Ident,    ThreadID, TaskData, IfVal,     lb,     ub,
+                     loadstep, NoGroup,  Sched,    GrainSize, TaskDup};
+
+    // taskloop runtime call
+    Function *TaskloopFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_taskloop);
+    Builder.CreateCall(TaskloopFn, Args);
+
+    // Emit the @__kmpc_end_taskgroup runtime call to end the taskgroup
+    Function *EndTaskgroupFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_end_taskgroup);
+    Builder.CreateCall(EndTaskgroupFn, {Ident, ThreadID});
+
+    StaleCI->eraseFromParent();
+
+    Builder.SetInsertPoint(TaskloopAllocaBB, TaskloopAllocaBB->begin());
+
+    if (HasShareds) {
+      LoadInst *Shareds = Builder.CreateLoad(VoidPtr, OutlinedFn.getArg(1));
+      OutlinedFn.getArg(1)->replaceUsesWithIf(
+          Shareds, [Shareds](Use &U) { return U.getUser() != Shareds; });
+    }
+
+    Value *IV = CLI->getIndVar();
+    Type *IVTy = IV->getType();
+    Constant *One = ConstantInt::get(IVTy, 1);
+
+    Value *task_lb = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
+                                             OutlinedFn.getArg(1), 5, "gep_lb");
+    Value *LowerBound = Builder.CreateLoad(IVTy, task_lb, "lb");
+
+    Value *task_ub = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
+                                             OutlinedFn.getArg(1), 6, "gep_ub");
+    Value *UpperBound = Builder.CreateLoad(IVTy, task_ub, "ub");
+
+    Builder.SetInsertPoint(CLI->getPreheader()->getTerminator());
+
+    Value *TripCountMinusOne = Builder.CreateSub(UpperBound, LowerBound);
+    Value *TripCount = Builder.CreateAdd(TripCountMinusOne, One, "trip_cnt");
+    // set the trip count in the CLI
+    CLI->setTripCount(TripCount);
+
+    Builder.SetInsertPoint(CLI->getBody(),
+                           CLI->getBody()->getFirstInsertionPt());
+
+    llvm::BasicBlock *Body = CLI->getBody();
+    for (llvm::Instruction &I : *Body) {
+      if (auto *Add = llvm::dyn_cast<llvm::BinaryOperator>(&I)) {
+        if (Add->getOpcode() == llvm::Instruction::Add) {
+          if (llvm::isa<llvm::BinaryOperator>(Add->getOperand(0))) {
+            // update the starting index of the loop
+            Add->setOperand(1, LowerBound);
+          }
+        }
+      }
+    }
+
+    for (Instruction *I : llvm::reverse(ToBeDeleted)) {
+      I->eraseFromParent();
+    }
+  };
+
+  addOutlineInfo(std::move(OI));
+  Builder.SetInsertPoint(TaskloopExitBB, TaskloopExitBB->begin());
+  return Builder.saveIP();
+}
+
 OpenMPIRBuilder::InsertPointOrErrorTy OpenMPIRBuilder::createTask(
     const LocationDescription &Loc, InsertPointTy AllocaIP,
     BodyGenCallbackTy BodyGenCB, bool Tied, Value *Final, Value *IfCondition,
diff --git a/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp b/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
index 8edec990eaaba..d69fcd3db0413 100644
--- a/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
+++ b/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
@@ -323,6 +323,18 @@ static LogicalResult checkImplementationStatus(Operation &op) {
     if (op.getDistScheduleChunkSize())
       result = todo("dist_schedule with chunk_size");
   };
+  auto checkFinal = [&todo](auto op, LogicalResult &result) {
+    if (op.getFinal())
+      result = todo("final");
+  };
+  auto checkGrainsize = [&todo](auto op, LogicalResult &result) {
+    if (op.getGrainsize())
+      result = todo("grainsize");
+  };
+  auto checkIf = [](auto op, LogicalResult &) {
+    if (op.getIfExpr())
+      op.emitWarning("if");
+  };
   auto checkHint = [](auto op, LogicalResult &) {
     if (op.getHint())
       op.emitWarning("hint clause discarded");
@@ -340,10 +352,22 @@ static LogicalResult checkImplementationStatus(Operation &op) {
     if (!op.getLinearVars().empty() || !op.getLinearStepVars().empty())
       result = todo("linear");
   };
+  auto checkMergeable = [&todo](auto op, LogicalResult &result) {
+    if (op.getMergeable())
+      result = todo("mergeable");
+  };
+  auto checkNogroup = [&todo](auto op, LogicalResult &result) {
+    if (op.getNogroup())
+      result = todo("nogroup");
+  };
   auto checkNowait = [&todo](auto op, LogicalResult &result) {
     if (op.getNowait())
       result = todo("nowait");
   };
+  auto checkNumTasks = [&todo](auto op, LogicalResult &result) {
+    if (op.getNumTasks())
+      result = todo("num_tasks");
+  };
   auto checkOrder = [&todo](auto op, LogicalResult &result) {
     if (op.getOrder() || op.getOrderMod())
       result = todo("order");
@@ -417,7 +441,15 @@ static LogicalResult checkImplementationStatus(Operation &op) {
         checkNowait(op, result);
       })
       .Case([&](omp::TaskloopOp op) {
-        // TODO: Add other clauses check
+        checkAllocate(op, result);
+        checkFinal(op, result);
+        checkGrainsize(op, result);
+        checkIf(op, result);
+        checkInReduction(op, result);
+        checkMergeable(op, result);
+        checkNogroup(op, result);
+        checkNumTasks(op, result);
+        checkReduction(op, result);
         checkUntied(op, result);
         checkPriority(op, result);
       })
@@ -2097,6 +2129,8 @@ class TaskContextStructManager {
   /// private decls.
   void createGEPsToPrivateVars();
 
+  llvm::Value *isAllocated();
+
   /// De-allocate the task context structure.
   void freeStructPtr();
 
@@ -2177,13 +2211,26 @@ void TaskContextStructManager::createGEPsToPrivateVars() {
   }
 }
 
+llvm::Value *TaskContextStructManager::isAllocated() {
+  if (!structPtr)
+    return nullptr;
+
+  return builder.CreateIsNotNull(structPtr);
+}
+
 void TaskContextStructManager::freeStructPtr() {
   if (!structPtr)
     return;
 
   llvm::IRBuilderBase::InsertPointGuard guard{builder};
-  // Ensure we don't put the call to free() after the terminator
-  builder.SetInsertPoint(builder.GetInsertBlock()->getTerminator());
+  llvm::BasicBlock *currentBlock = builder.GetInsertBlock();
+  if (currentBlock->getTerminator()) {
+    // Ensure we don't put the call to free() after the terminator
+    builder.SetInsertPoint(currentBlock->getTerminator());
+  } else {
+    // Insert the call to free() at the end of the current block
+    builder.SetInsertPoint(currentBlock);
+  }
   builder.CreateFree(structPtr);
 }
 
@@ -2419,6 +2466,207 @@ convertOmpTaskOp(omp::TaskOp taskOp, llvm::IRBuilderBase &builder,
   return success();
 }
 
+// Converts an OpenMP taskloop construct into LLVM IR using OpenMPIRBuilder.
+static LogicalResult
+convertOmpTaskloopOp(Operation &opInst, llvm::IRBuilderBase &builder,
+                     LLVM::ModuleTranslation &moduleTranslation) {
+  using InsertPointTy = llvm::OpenMPIRBuilder::InsertPointTy;
+  auto taskloopOp = cast<omp::TaskloopOp>(opInst);
+  if (failed(checkImplementationStatus(opInst)))
+    return failure();
+
+  // It stores the pointer of allocated firstprivate copies,
+  // which can be used later for freeing the allocated space.
+  SmallVector<llvm::Value *> llvmFirstPrivateVars;
+  PrivateVarsInfo privateVarsInfo(taskloopOp);
+  TaskContextStructManager taskStructMgr{builder, moduleTranslation,
+                                         privateVarsInfo.privatizers};
+
+  llvm::OpenMPIRBuilder::InsertPointTy allocaIP =
+      findAllocaInsertPoint(builder, moduleTranslation);
+
+  assert(builder.GetInsertPoint() == builder.GetInsertBlock()->end());
+  llvm::BasicBlock *taskloopStartBlock = llvm::BasicBlock::Create(
+      builder.getContext(), "omp.taskloop.start",
+      /*Parent=*/builder.GetInsertBlock()->getParent());
+  llvm::Instruction *branchToTaskloopStartBlock =
+      builder.CreateBr(taskloopStartBlock);
+  builder.SetInsertPoint(branchToTaskloopStartBlock);
+
+  llvm::BasicBlock *copyBlock =
+      splitBB(builder, /*CreateBranch=*/true, "omp.private.copy");
+  llvm::BasicBlock *initBlock =
+      splitBB(builder, /*CreateBranch=*/true, "omp.private.init");
+
+  LLVM::ModuleTranslation::SaveStack<OpenMPAllocaStackFrame> frame(
+      moduleTranslation, allocaIP);
+
+  // Allocate and initialize private variables
+  builder.SetInsertPoint(initBlock->getTerminator());
+
+  taskStructMgr.generateTaskContextStruct();
+  taskStructMgr.createGEPsToPrivateVars();
+
+  llvmFirstPrivateVars.resize(privateVarsInfo.blockArgs.size());
+  int index = 0;
+
+  for (auto [privDecl, mlirPrivVar, blockArg, llvmPrivateVarAlloc] :
+       llvm::zip_equal(privateVarsInfo.privatizers, privateVarsInfo.mlirVars,
+                       privateVarsInfo.blockArgs,
+                       taskStructMgr.getLLVMPrivateVarGEPs())) {
+    // To be handled inside the taskloop.
+    if (!privDecl.readsFromMold())
+      continue;
+    assert(llvmPrivateVarAlloc &&
+           "reads from mold so shouldn't have been skipped");
+
+    llvm::Expected<llvm::Value *> privateVarOrErr =
+        initPrivateVar(builder, moduleTranslation, privDecl, mlirPrivVar,
+                       blockArg, llvmPrivateVarAlloc, initBlock);
+    if (!privateVarOrErr)
+      return handleError(privateVarOrErr, *taskloopOp.getOperation());
+
+    llvmFirstPrivateVars[index++] = privateVarOrErr.get();
+
+    llvm::IRBuilderBase::InsertPointGuard guard(builder);
+    builder.SetInsertPoint(builder.GetInsertBlock()->getTerminator());
+
+    if ((privateVarOrErr.get() != llvmPrivateVarAlloc) &&
+        !mlir::isa<LLVM::LLVMPointerType>(blockArg.getType())) {
+      builder.CreateStore(privateVarOrErr.get(), llvmPrivateVarAlloc);
+      // Load it so we have the value pointed to by the GEP
+      llvmPrivateVarAlloc = builder.CreateLoad(privateVarOrErr.get()->getType(),
+                                               llvmPrivateVarAlloc);
+    }
+    assert(llvmPrivateVarAlloc->getType() ==
+           moduleTranslation.convertType(blockArg.getType()));
+  }
+
+  // firstprivate copy region
+  setInsertPointForPossiblyEmptyBlock(builder, copyBlock);
+  if (failed(copyFirstPrivateVars(
+          taskloopOp, builder, moduleTranslation, privateVarsInfo.mlirVars,
+          taskStructMgr.getLLVMPrivateVarGEPs(), privateVarsInfo.privatizers,
+          taskloopOp.getPrivateNeedsBarrier())))
+    return llvm::failure();
+
+  // Set up inserttion point for call to createTaskloop()
+  builder.SetInsertPoint(taskloopStartBlock);
+
+  auto bodyCB = [&](InsertPointTy allocaIP,
+                    InsertPointTy codegenIP) -> llvm::Error {
+    // Save the alloca insertion point on ModuleTranslation stack for use in
+    // nested regions.
+    LLVM::ModuleTranslation::SaveStack<OpenMPAllocaStackFrame> frame(
+        moduleTranslation, allocaIP);
+
+    // translate the body of the taskloop:
+    builder.restoreIP(codegenIP);
+
+    llvm::BasicBlock *privInitBlock = nullptr;
+    privateVarsInfo.llvmVars.resize(privateVarsInfo.blockArgs.size());
+    for (auto [i, zip] : llvm::enumerate(llvm::zip_equal(
+             privateVarsInfo.blockArgs, privateVarsInfo.privatizers,
+             privateVarsInfo.mlirVars))) {
+      auto [blockArg, privDecl, mlirPrivVar] = zip;
+      // This is handled before the task executes
+      if (privDecl.readsFromMold())
+        continue;
+
+      llvm::IRBuilderBase::InsertPointGuard guard(builder);
+      llvm::Type *llvmAllocType =
+          moduleTranslation.convertType(privDecl.getType());
+      builder.SetInsertPoint(allocaIP.getBlock()->getTerminator());
+      llvm::Value *llvmPrivateVar = builder.CreateAlloca(
+          llvmAllocType, /*ArraySize=*/nullptr, "omp.private.alloc");
+
+      llvm::Expected<llvm::Value *> privateVarOrError =
+          initPrivateVar(builder, moduleTranslation, privDecl, mlirPrivVar,
+                         blockArg, llvmPrivateVar, privInitBlock);
+      if (!privateVarOrError)
+        return privateVarOrError.takeError();
+      moduleTranslation.mapValue(blockArg, privateVarOrError.get());
+      privateVarsInfo.llvmVars[i] = privateVarOrError.get();
+      // Add private var to  llvmFirstPrivateVars
+      llvmFirstPrivateVars[index++] = privateVarOrError.get();
+    }
+
+    taskStructMgr.createGEPsToPrivateVars();
+    for (auto [i, llvmPrivVar] :
+         llvm::enumerate(taskStructMgr.getLLVMPrivateVarGEPs())) {
+      if (!llvmPrivVar) {
...
[truncated]

@llvmbot
Copy link
Member

llvmbot commented Nov 7, 2025

@llvm/pr-subscribers-flang-openmp

Author: Kaviya Rajendiran (kaviya2510)

Changes

Added translation support for taskloop construct.


Patch is 32.13 KiB, truncated to 20.00 KiB below, full version: https://github.com/llvm/llvm-project/pull/166903.diff

6 Files Affected:

  • (modified) llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h (+16)
  • (modified) llvm/include/llvm/Frontend/OpenMP/OMPKinds.def (+1)
  • (modified) llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp (+199)
  • (modified) mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp (+254-3)
  • (added) mlir/test/Target/LLVMIR/openmp-taskloop.mlir (+151)
  • (modified) mlir/test/Target/LLVMIR/openmp-todo.mlir (+1-14)
diff --git a/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h b/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
index b3d7ab4acf303..18828380abd32 100644
--- a/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
+++ b/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
@@ -1359,6 +1359,22 @@ class OpenMPIRBuilder {
         : DepKind(DepKind), DepValueType(DepValueType), DepVal(DepVal) {}
   };
 
+  /// Generator for `#omp taskloop`
+  ///
+  /// \param Loc The location where the taskloop construct was encountered.
+  /// \param AllocaIP The insertion point to be used for alloca instructions.
+  /// \param BodyGenCB Callback that will generate the region code.
+  /// \param LoopInfo Callback that return the CLI
+  /// \param LBVal Lowerbound value of loop
+  /// \param UBVal Upperbound value of loop
+  /// \param StepVal Step value of loop
+  /// \param Tied True if the task is tied, false if the task is untied.
+  LLVM_ABI InsertPointOrErrorTy createTaskloop(
+      const LocationDescription &Loc, InsertPointTy AllocaIP,
+      BodyGenCallbackTy BodyGenCB,
+      llvm::function_ref<llvm::Expected<llvm::CanonicalLoopInfo *>()> LoopInfo,
+      Value *LBVal, Value *UBVal, Value *StepVal, bool Tied = true);
+
   /// Generator for `#omp task`
   ///
   /// \param Loc The location where the task construct was encountered.
diff --git a/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def b/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
index 46b3d53a4b408..032495dfe9d61 100644
--- a/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
+++ b/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
@@ -95,6 +95,7 @@ __OMP_STRUCT_TYPE(KernelArgs, __tgt_kernel_arguments, false, Int32, Int32, VoidP
 __OMP_STRUCT_TYPE(AsyncInfo, __tgt_async_info, false, Int8Ptr)
 __OMP_STRUCT_TYPE(DependInfo, kmp_dep_info, false, SizeTy, SizeTy, Int8)
 __OMP_STRUCT_TYPE(Task, kmp_task_ompbuilder_t, false, VoidPtr, VoidPtr, Int32, VoidPtr, VoidPtr)
+__OMP_STRUCT_TYPE(Taskloop, kmp_task_info, false, VoidPtr, VoidPtr, Int32, VoidPtr, VoidPtr, Int64, Int64, Int64)
 __OMP_STRUCT_TYPE(ConfigurationEnvironment, ConfigurationEnvironmentTy, false,
                   Int8, Int8, Int8, Int32, Int32, Int32, Int32, Int32, Int32)
 __OMP_STRUCT_TYPE(DynamicEnvironment, DynamicEnvironmentTy, false, Int16)
diff --git a/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp b/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
index fff9a815e5368..e88e722b1370e 100644
--- a/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
+++ b/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
@@ -1933,6 +1933,205 @@ static Value *emitTaskDependencies(
   return DepArray;
 }
 
+OpenMPIRBuilder::InsertPointOrErrorTy OpenMPIRBuilder::createTaskloop(
+    const LocationDescription &Loc, InsertPointTy AllocaIP,
+    BodyGenCallbackTy BodyGenCB,
+    llvm::function_ref<llvm::Expected<llvm::CanonicalLoopInfo *>()> loopInfo,
+    Value *LBVal, Value *UBVal, Value *StepVal, bool Tied) {
+
+  if (!updateToLocation(Loc))
+    return InsertPointTy();
+
+  uint32_t SrcLocStrSize;
+  Constant *SrcLocStr = getOrCreateSrcLocStr(Loc, SrcLocStrSize);
+  Value *Ident = getOrCreateIdent(SrcLocStr, SrcLocStrSize);
+
+  BasicBlock *TaskloopExitBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.exit");
+  BasicBlock *TaskloopBodyBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.body");
+  BasicBlock *TaskloopAllocaBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.alloca");
+
+  InsertPointTy TaskloopAllocaIP =
+      InsertPointTy(TaskloopAllocaBB, TaskloopAllocaBB->begin());
+  InsertPointTy TaskloopBodyIP =
+      InsertPointTy(TaskloopBodyBB, TaskloopBodyBB->begin());
+
+  if (Error Err = BodyGenCB(TaskloopAllocaIP, TaskloopBodyIP))
+    return Err;
+
+  llvm::Expected<llvm::CanonicalLoopInfo *> result = loopInfo();
+  if (!result) {
+    return result.takeError();
+  }
+
+  llvm::CanonicalLoopInfo *CLI = result.get();
+  OutlineInfo OI;
+  OI.EntryBB = TaskloopAllocaBB;
+  OI.OuterAllocaBB = AllocaIP.getBlock();
+  OI.ExitBB = TaskloopExitBB;
+
+  // Add the thread ID argument.
+  SmallVector<Instruction *, 4> ToBeDeleted;
+  // dummy instruction to be used as a fake argument
+  OI.ExcludeArgsFromAggregate.push_back(createFakeIntVal(
+      Builder, AllocaIP, ToBeDeleted, TaskloopAllocaIP, "global.tid", false));
+
+  OI.PostOutlineCB = [this, Ident, LBVal, UBVal, StepVal, Tied,
+                      TaskloopAllocaBB, CLI, Loc,
+                      ToBeDeleted](Function &OutlinedFn) mutable {
+    // Replace the Stale CI by appropriate RTL function call.
+    assert(OutlinedFn.hasOneUse() &&
+           "there must be a single user for the outlined function");
+    CallInst *StaleCI = cast<CallInst>(OutlinedFn.user_back());
+
+    // HasShareds is true if any variables are captured in the outlined region,
+    // false otherwise.
+    bool HasShareds = StaleCI->arg_size() > 1;
+    Builder.SetInsertPoint(StaleCI);
+
+    // Gather the arguments for emitting the runtime call for
+    // @__kmpc_omp_task_alloc
+    Function *TaskAllocFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_omp_task_alloc);
+
+    Value *ThreadID = getOrCreateThreadID(Ident);
+
+    // Emit runtime call for @__kmpc_taskgroup
+    Function *TaskgroupFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_taskgroup);
+    Builder.CreateCall(TaskgroupFn, {Ident, ThreadID});
+
+    // The flags are set to 1 if the task is tied, 0 otherwise.
+    Value *Flags = Builder.getInt32(Tied);
+
+    Value *TaskSize = Builder.getInt64(
+        divideCeil(M.getDataLayout().getTypeSizeInBits(Taskloop), 8));
+
+    Value *SharedsSize = Builder.getInt64(0);
+    if (HasShareds) {
+      AllocaInst *ArgStructAlloca =
+          dyn_cast<AllocaInst>(StaleCI->getArgOperand(1));
+      assert(ArgStructAlloca &&
+             "Unable to find the alloca instruction corresponding to arguments "
+             "for extracted function");
+      StructType *ArgStructType =
+          dyn_cast<StructType>(ArgStructAlloca->getAllocatedType());
+      assert(ArgStructType && "Unable to find struct type corresponding to "
+                              "arguments for extracted function");
+      SharedsSize =
+          Builder.getInt64(M.getDataLayout().getTypeStoreSize(ArgStructType));
+    }
+
+    // Emit the @__kmpc_omp_task_alloc runtime call
+    // The runtime call returns a pointer to an area where the task captured
+    // variables must be copied before the task is run (TaskData)
+    CallInst *TaskData = Builder.CreateCall(
+        TaskAllocFn, {/*loc_ref=*/Ident, /*gtid=*/ThreadID, /*flags=*/Flags,
+                      /*sizeof_task=*/TaskSize, /*sizeof_shared=*/SharedsSize,
+                      /*task_func=*/&OutlinedFn});
+
+    // Get the pointer to loop lb, ub, step from task ptr
+    // and set up the lowerbound,upperbound and step values
+    llvm::Value *lb =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 5);
+    //  Value *LbVal_ext = Builder.CreateSExt(LBVal, Builder.getInt64Ty());
+    Builder.CreateStore(LBVal, lb);
+
+    llvm::Value *ub =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 6);
+    Builder.CreateStore(UBVal, ub);
+
+    llvm::Value *step =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 7);
+    Value *Step_ext = Builder.CreateSExt(StepVal, Builder.getInt64Ty());
+    Builder.CreateStore(Step_ext, step);
+    llvm::Value *loadstep = Builder.CreateLoad(Builder.getInt64Ty(), step);
+
+    if (HasShareds) {
+      Value *Shareds = StaleCI->getArgOperand(1);
+      Align Alignment = TaskData->getPointerAlignment(M.getDataLayout());
+      Value *TaskShareds = Builder.CreateLoad(VoidPtr, TaskData);
+      Builder.CreateMemCpy(TaskShareds, Alignment, Shareds, Alignment,
+                           SharedsSize);
+    }
+
+    // set up the arguments for emitting kmpc_taskloop runtime call
+    // setting default values for ifval, nogroup, sched, grainsize, task_dup
+    Value *IfVal = Builder.getInt32(1);
+    Value *NoGroup = Builder.getInt32(1);
+    Value *Sched = Builder.getInt32(0);
+    Value *GrainSize = Builder.getInt64(0);
+    Value *TaskDup = Constant::getNullValue(Builder.getPtrTy());
+
+    Value *Args[] = {Ident,    ThreadID, TaskData, IfVal,     lb,     ub,
+                     loadstep, NoGroup,  Sched,    GrainSize, TaskDup};
+
+    // taskloop runtime call
+    Function *TaskloopFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_taskloop);
+    Builder.CreateCall(TaskloopFn, Args);
+
+    // Emit the @__kmpc_end_taskgroup runtime call to end the taskgroup
+    Function *EndTaskgroupFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_end_taskgroup);
+    Builder.CreateCall(EndTaskgroupFn, {Ident, ThreadID});
+
+    StaleCI->eraseFromParent();
+
+    Builder.SetInsertPoint(TaskloopAllocaBB, TaskloopAllocaBB->begin());
+
+    if (HasShareds) {
+      LoadInst *Shareds = Builder.CreateLoad(VoidPtr, OutlinedFn.getArg(1));
+      OutlinedFn.getArg(1)->replaceUsesWithIf(
+          Shareds, [Shareds](Use &U) { return U.getUser() != Shareds; });
+    }
+
+    Value *IV = CLI->getIndVar();
+    Type *IVTy = IV->getType();
+    Constant *One = ConstantInt::get(IVTy, 1);
+
+    Value *task_lb = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
+                                             OutlinedFn.getArg(1), 5, "gep_lb");
+    Value *LowerBound = Builder.CreateLoad(IVTy, task_lb, "lb");
+
+    Value *task_ub = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
+                                             OutlinedFn.getArg(1), 6, "gep_ub");
+    Value *UpperBound = Builder.CreateLoad(IVTy, task_ub, "ub");
+
+    Builder.SetInsertPoint(CLI->getPreheader()->getTerminator());
+
+    Value *TripCountMinusOne = Builder.CreateSub(UpperBound, LowerBound);
+    Value *TripCount = Builder.CreateAdd(TripCountMinusOne, One, "trip_cnt");
+    // set the trip count in the CLI
+    CLI->setTripCount(TripCount);
+
+    Builder.SetInsertPoint(CLI->getBody(),
+                           CLI->getBody()->getFirstInsertionPt());
+
+    llvm::BasicBlock *Body = CLI->getBody();
+    for (llvm::Instruction &I : *Body) {
+      if (auto *Add = llvm::dyn_cast<llvm::BinaryOperator>(&I)) {
+        if (Add->getOpcode() == llvm::Instruction::Add) {
+          if (llvm::isa<llvm::BinaryOperator>(Add->getOperand(0))) {
+            // update the starting index of the loop
+            Add->setOperand(1, LowerBound);
+          }
+        }
+      }
+    }
+
+    for (Instruction *I : llvm::reverse(ToBeDeleted)) {
+      I->eraseFromParent();
+    }
+  };
+
+  addOutlineInfo(std::move(OI));
+  Builder.SetInsertPoint(TaskloopExitBB, TaskloopExitBB->begin());
+  return Builder.saveIP();
+}
+
 OpenMPIRBuilder::InsertPointOrErrorTy OpenMPIRBuilder::createTask(
     const LocationDescription &Loc, InsertPointTy AllocaIP,
     BodyGenCallbackTy BodyGenCB, bool Tied, Value *Final, Value *IfCondition,
diff --git a/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp b/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
index 8edec990eaaba..d69fcd3db0413 100644
--- a/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
+++ b/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
@@ -323,6 +323,18 @@ static LogicalResult checkImplementationStatus(Operation &op) {
     if (op.getDistScheduleChunkSize())
       result = todo("dist_schedule with chunk_size");
   };
+  auto checkFinal = [&todo](auto op, LogicalResult &result) {
+    if (op.getFinal())
+      result = todo("final");
+  };
+  auto checkGrainsize = [&todo](auto op, LogicalResult &result) {
+    if (op.getGrainsize())
+      result = todo("grainsize");
+  };
+  auto checkIf = [](auto op, LogicalResult &) {
+    if (op.getIfExpr())
+      op.emitWarning("if");
+  };
   auto checkHint = [](auto op, LogicalResult &) {
     if (op.getHint())
       op.emitWarning("hint clause discarded");
@@ -340,10 +352,22 @@ static LogicalResult checkImplementationStatus(Operation &op) {
     if (!op.getLinearVars().empty() || !op.getLinearStepVars().empty())
       result = todo("linear");
   };
+  auto checkMergeable = [&todo](auto op, LogicalResult &result) {
+    if (op.getMergeable())
+      result = todo("mergeable");
+  };
+  auto checkNogroup = [&todo](auto op, LogicalResult &result) {
+    if (op.getNogroup())
+      result = todo("nogroup");
+  };
   auto checkNowait = [&todo](auto op, LogicalResult &result) {
     if (op.getNowait())
       result = todo("nowait");
   };
+  auto checkNumTasks = [&todo](auto op, LogicalResult &result) {
+    if (op.getNumTasks())
+      result = todo("num_tasks");
+  };
   auto checkOrder = [&todo](auto op, LogicalResult &result) {
     if (op.getOrder() || op.getOrderMod())
       result = todo("order");
@@ -417,7 +441,15 @@ static LogicalResult checkImplementationStatus(Operation &op) {
         checkNowait(op, result);
       })
       .Case([&](omp::TaskloopOp op) {
-        // TODO: Add other clauses check
+        checkAllocate(op, result);
+        checkFinal(op, result);
+        checkGrainsize(op, result);
+        checkIf(op, result);
+        checkInReduction(op, result);
+        checkMergeable(op, result);
+        checkNogroup(op, result);
+        checkNumTasks(op, result);
+        checkReduction(op, result);
         checkUntied(op, result);
         checkPriority(op, result);
       })
@@ -2097,6 +2129,8 @@ class TaskContextStructManager {
   /// private decls.
   void createGEPsToPrivateVars();
 
+  llvm::Value *isAllocated();
+
   /// De-allocate the task context structure.
   void freeStructPtr();
 
@@ -2177,13 +2211,26 @@ void TaskContextStructManager::createGEPsToPrivateVars() {
   }
 }
 
+llvm::Value *TaskContextStructManager::isAllocated() {
+  if (!structPtr)
+    return nullptr;
+
+  return builder.CreateIsNotNull(structPtr);
+}
+
 void TaskContextStructManager::freeStructPtr() {
   if (!structPtr)
     return;
 
   llvm::IRBuilderBase::InsertPointGuard guard{builder};
-  // Ensure we don't put the call to free() after the terminator
-  builder.SetInsertPoint(builder.GetInsertBlock()->getTerminator());
+  llvm::BasicBlock *currentBlock = builder.GetInsertBlock();
+  if (currentBlock->getTerminator()) {
+    // Ensure we don't put the call to free() after the terminator
+    builder.SetInsertPoint(currentBlock->getTerminator());
+  } else {
+    // Insert the call to free() at the end of the current block
+    builder.SetInsertPoint(currentBlock);
+  }
   builder.CreateFree(structPtr);
 }
 
@@ -2419,6 +2466,207 @@ convertOmpTaskOp(omp::TaskOp taskOp, llvm::IRBuilderBase &builder,
   return success();
 }
 
+// Converts an OpenMP taskloop construct into LLVM IR using OpenMPIRBuilder.
+static LogicalResult
+convertOmpTaskloopOp(Operation &opInst, llvm::IRBuilderBase &builder,
+                     LLVM::ModuleTranslation &moduleTranslation) {
+  using InsertPointTy = llvm::OpenMPIRBuilder::InsertPointTy;
+  auto taskloopOp = cast<omp::TaskloopOp>(opInst);
+  if (failed(checkImplementationStatus(opInst)))
+    return failure();
+
+  // It stores the pointer of allocated firstprivate copies,
+  // which can be used later for freeing the allocated space.
+  SmallVector<llvm::Value *> llvmFirstPrivateVars;
+  PrivateVarsInfo privateVarsInfo(taskloopOp);
+  TaskContextStructManager taskStructMgr{builder, moduleTranslation,
+                                         privateVarsInfo.privatizers};
+
+  llvm::OpenMPIRBuilder::InsertPointTy allocaIP =
+      findAllocaInsertPoint(builder, moduleTranslation);
+
+  assert(builder.GetInsertPoint() == builder.GetInsertBlock()->end());
+  llvm::BasicBlock *taskloopStartBlock = llvm::BasicBlock::Create(
+      builder.getContext(), "omp.taskloop.start",
+      /*Parent=*/builder.GetInsertBlock()->getParent());
+  llvm::Instruction *branchToTaskloopStartBlock =
+      builder.CreateBr(taskloopStartBlock);
+  builder.SetInsertPoint(branchToTaskloopStartBlock);
+
+  llvm::BasicBlock *copyBlock =
+      splitBB(builder, /*CreateBranch=*/true, "omp.private.copy");
+  llvm::BasicBlock *initBlock =
+      splitBB(builder, /*CreateBranch=*/true, "omp.private.init");
+
+  LLVM::ModuleTranslation::SaveStack<OpenMPAllocaStackFrame> frame(
+      moduleTranslation, allocaIP);
+
+  // Allocate and initialize private variables
+  builder.SetInsertPoint(initBlock->getTerminator());
+
+  taskStructMgr.generateTaskContextStruct();
+  taskStructMgr.createGEPsToPrivateVars();
+
+  llvmFirstPrivateVars.resize(privateVarsInfo.blockArgs.size());
+  int index = 0;
+
+  for (auto [privDecl, mlirPrivVar, blockArg, llvmPrivateVarAlloc] :
+       llvm::zip_equal(privateVarsInfo.privatizers, privateVarsInfo.mlirVars,
+                       privateVarsInfo.blockArgs,
+                       taskStructMgr.getLLVMPrivateVarGEPs())) {
+    // To be handled inside the taskloop.
+    if (!privDecl.readsFromMold())
+      continue;
+    assert(llvmPrivateVarAlloc &&
+           "reads from mold so shouldn't have been skipped");
+
+    llvm::Expected<llvm::Value *> privateVarOrErr =
+        initPrivateVar(builder, moduleTranslation, privDecl, mlirPrivVar,
+                       blockArg, llvmPrivateVarAlloc, initBlock);
+    if (!privateVarOrErr)
+      return handleError(privateVarOrErr, *taskloopOp.getOperation());
+
+    llvmFirstPrivateVars[index++] = privateVarOrErr.get();
+
+    llvm::IRBuilderBase::InsertPointGuard guard(builder);
+    builder.SetInsertPoint(builder.GetInsertBlock()->getTerminator());
+
+    if ((privateVarOrErr.get() != llvmPrivateVarAlloc) &&
+        !mlir::isa<LLVM::LLVMPointerType>(blockArg.getType())) {
+      builder.CreateStore(privateVarOrErr.get(), llvmPrivateVarAlloc);
+      // Load it so we have the value pointed to by the GEP
+      llvmPrivateVarAlloc = builder.CreateLoad(privateVarOrErr.get()->getType(),
+                                               llvmPrivateVarAlloc);
+    }
+    assert(llvmPrivateVarAlloc->getType() ==
+           moduleTranslation.convertType(blockArg.getType()));
+  }
+
+  // firstprivate copy region
+  setInsertPointForPossiblyEmptyBlock(builder, copyBlock);
+  if (failed(copyFirstPrivateVars(
+          taskloopOp, builder, moduleTranslation, privateVarsInfo.mlirVars,
+          taskStructMgr.getLLVMPrivateVarGEPs(), privateVarsInfo.privatizers,
+          taskloopOp.getPrivateNeedsBarrier())))
+    return llvm::failure();
+
+  // Set up inserttion point for call to createTaskloop()
+  builder.SetInsertPoint(taskloopStartBlock);
+
+  auto bodyCB = [&](InsertPointTy allocaIP,
+                    InsertPointTy codegenIP) -> llvm::Error {
+    // Save the alloca insertion point on ModuleTranslation stack for use in
+    // nested regions.
+    LLVM::ModuleTranslation::SaveStack<OpenMPAllocaStackFrame> frame(
+        moduleTranslation, allocaIP);
+
+    // translate the body of the taskloop:
+    builder.restoreIP(codegenIP);
+
+    llvm::BasicBlock *privInitBlock = nullptr;
+    privateVarsInfo.llvmVars.resize(privateVarsInfo.blockArgs.size());
+    for (auto [i, zip] : llvm::enumerate(llvm::zip_equal(
+             privateVarsInfo.blockArgs, privateVarsInfo.privatizers,
+             privateVarsInfo.mlirVars))) {
+      auto [blockArg, privDecl, mlirPrivVar] = zip;
+      // This is handled before the task executes
+      if (privDecl.readsFromMold())
+        continue;
+
+      llvm::IRBuilderBase::InsertPointGuard guard(builder);
+      llvm::Type *llvmAllocType =
+          moduleTranslation.convertType(privDecl.getType());
+      builder.SetInsertPoint(allocaIP.getBlock()->getTerminator());
+      llvm::Value *llvmPrivateVar = builder.CreateAlloca(
+          llvmAllocType, /*ArraySize=*/nullptr, "omp.private.alloc");
+
+      llvm::Expected<llvm::Value *> privateVarOrError =
+          initPrivateVar(builder, moduleTranslation, privDecl, mlirPrivVar,
+                         blockArg, llvmPrivateVar, privInitBlock);
+      if (!privateVarOrError)
+        return privateVarOrError.takeError();
+      moduleTranslation.mapValue(blockArg, privateVarOrError.get());
+      privateVarsInfo.llvmVars[i] = privateVarOrError.get();
+      // Add private var to  llvmFirstPrivateVars
+      llvmFirstPrivateVars[index++] = privateVarOrError.get();
+    }
+
+    taskStructMgr.createGEPsToPrivateVars();
+    for (auto [i, llvmPrivVar] :
+         llvm::enumerate(taskStructMgr.getLLVMPrivateVarGEPs())) {
+      if (!llvmPrivVar) {
...
[truncated]

@llvmbot
Copy link
Member

llvmbot commented Nov 7, 2025

@llvm/pr-subscribers-mlir

Author: Kaviya Rajendiran (kaviya2510)

Changes

Added translation support for taskloop construct.


Patch is 32.13 KiB, truncated to 20.00 KiB below, full version: https://github.com/llvm/llvm-project/pull/166903.diff

6 Files Affected:

  • (modified) llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h (+16)
  • (modified) llvm/include/llvm/Frontend/OpenMP/OMPKinds.def (+1)
  • (modified) llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp (+199)
  • (modified) mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp (+254-3)
  • (added) mlir/test/Target/LLVMIR/openmp-taskloop.mlir (+151)
  • (modified) mlir/test/Target/LLVMIR/openmp-todo.mlir (+1-14)
diff --git a/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h b/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
index b3d7ab4acf303..18828380abd32 100644
--- a/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
+++ b/llvm/include/llvm/Frontend/OpenMP/OMPIRBuilder.h
@@ -1359,6 +1359,22 @@ class OpenMPIRBuilder {
         : DepKind(DepKind), DepValueType(DepValueType), DepVal(DepVal) {}
   };
 
+  /// Generator for `#omp taskloop`
+  ///
+  /// \param Loc The location where the taskloop construct was encountered.
+  /// \param AllocaIP The insertion point to be used for alloca instructions.
+  /// \param BodyGenCB Callback that will generate the region code.
+  /// \param LoopInfo Callback that return the CLI
+  /// \param LBVal Lowerbound value of loop
+  /// \param UBVal Upperbound value of loop
+  /// \param StepVal Step value of loop
+  /// \param Tied True if the task is tied, false if the task is untied.
+  LLVM_ABI InsertPointOrErrorTy createTaskloop(
+      const LocationDescription &Loc, InsertPointTy AllocaIP,
+      BodyGenCallbackTy BodyGenCB,
+      llvm::function_ref<llvm::Expected<llvm::CanonicalLoopInfo *>()> LoopInfo,
+      Value *LBVal, Value *UBVal, Value *StepVal, bool Tied = true);
+
   /// Generator for `#omp task`
   ///
   /// \param Loc The location where the task construct was encountered.
diff --git a/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def b/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
index 46b3d53a4b408..032495dfe9d61 100644
--- a/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
+++ b/llvm/include/llvm/Frontend/OpenMP/OMPKinds.def
@@ -95,6 +95,7 @@ __OMP_STRUCT_TYPE(KernelArgs, __tgt_kernel_arguments, false, Int32, Int32, VoidP
 __OMP_STRUCT_TYPE(AsyncInfo, __tgt_async_info, false, Int8Ptr)
 __OMP_STRUCT_TYPE(DependInfo, kmp_dep_info, false, SizeTy, SizeTy, Int8)
 __OMP_STRUCT_TYPE(Task, kmp_task_ompbuilder_t, false, VoidPtr, VoidPtr, Int32, VoidPtr, VoidPtr)
+__OMP_STRUCT_TYPE(Taskloop, kmp_task_info, false, VoidPtr, VoidPtr, Int32, VoidPtr, VoidPtr, Int64, Int64, Int64)
 __OMP_STRUCT_TYPE(ConfigurationEnvironment, ConfigurationEnvironmentTy, false,
                   Int8, Int8, Int8, Int32, Int32, Int32, Int32, Int32, Int32)
 __OMP_STRUCT_TYPE(DynamicEnvironment, DynamicEnvironmentTy, false, Int16)
diff --git a/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp b/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
index fff9a815e5368..e88e722b1370e 100644
--- a/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
+++ b/llvm/lib/Frontend/OpenMP/OMPIRBuilder.cpp
@@ -1933,6 +1933,205 @@ static Value *emitTaskDependencies(
   return DepArray;
 }
 
+OpenMPIRBuilder::InsertPointOrErrorTy OpenMPIRBuilder::createTaskloop(
+    const LocationDescription &Loc, InsertPointTy AllocaIP,
+    BodyGenCallbackTy BodyGenCB,
+    llvm::function_ref<llvm::Expected<llvm::CanonicalLoopInfo *>()> loopInfo,
+    Value *LBVal, Value *UBVal, Value *StepVal, bool Tied) {
+
+  if (!updateToLocation(Loc))
+    return InsertPointTy();
+
+  uint32_t SrcLocStrSize;
+  Constant *SrcLocStr = getOrCreateSrcLocStr(Loc, SrcLocStrSize);
+  Value *Ident = getOrCreateIdent(SrcLocStr, SrcLocStrSize);
+
+  BasicBlock *TaskloopExitBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.exit");
+  BasicBlock *TaskloopBodyBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.body");
+  BasicBlock *TaskloopAllocaBB =
+      splitBB(Builder, /*CreateBranch=*/true, "taskloop.alloca");
+
+  InsertPointTy TaskloopAllocaIP =
+      InsertPointTy(TaskloopAllocaBB, TaskloopAllocaBB->begin());
+  InsertPointTy TaskloopBodyIP =
+      InsertPointTy(TaskloopBodyBB, TaskloopBodyBB->begin());
+
+  if (Error Err = BodyGenCB(TaskloopAllocaIP, TaskloopBodyIP))
+    return Err;
+
+  llvm::Expected<llvm::CanonicalLoopInfo *> result = loopInfo();
+  if (!result) {
+    return result.takeError();
+  }
+
+  llvm::CanonicalLoopInfo *CLI = result.get();
+  OutlineInfo OI;
+  OI.EntryBB = TaskloopAllocaBB;
+  OI.OuterAllocaBB = AllocaIP.getBlock();
+  OI.ExitBB = TaskloopExitBB;
+
+  // Add the thread ID argument.
+  SmallVector<Instruction *, 4> ToBeDeleted;
+  // dummy instruction to be used as a fake argument
+  OI.ExcludeArgsFromAggregate.push_back(createFakeIntVal(
+      Builder, AllocaIP, ToBeDeleted, TaskloopAllocaIP, "global.tid", false));
+
+  OI.PostOutlineCB = [this, Ident, LBVal, UBVal, StepVal, Tied,
+                      TaskloopAllocaBB, CLI, Loc,
+                      ToBeDeleted](Function &OutlinedFn) mutable {
+    // Replace the Stale CI by appropriate RTL function call.
+    assert(OutlinedFn.hasOneUse() &&
+           "there must be a single user for the outlined function");
+    CallInst *StaleCI = cast<CallInst>(OutlinedFn.user_back());
+
+    // HasShareds is true if any variables are captured in the outlined region,
+    // false otherwise.
+    bool HasShareds = StaleCI->arg_size() > 1;
+    Builder.SetInsertPoint(StaleCI);
+
+    // Gather the arguments for emitting the runtime call for
+    // @__kmpc_omp_task_alloc
+    Function *TaskAllocFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_omp_task_alloc);
+
+    Value *ThreadID = getOrCreateThreadID(Ident);
+
+    // Emit runtime call for @__kmpc_taskgroup
+    Function *TaskgroupFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_taskgroup);
+    Builder.CreateCall(TaskgroupFn, {Ident, ThreadID});
+
+    // The flags are set to 1 if the task is tied, 0 otherwise.
+    Value *Flags = Builder.getInt32(Tied);
+
+    Value *TaskSize = Builder.getInt64(
+        divideCeil(M.getDataLayout().getTypeSizeInBits(Taskloop), 8));
+
+    Value *SharedsSize = Builder.getInt64(0);
+    if (HasShareds) {
+      AllocaInst *ArgStructAlloca =
+          dyn_cast<AllocaInst>(StaleCI->getArgOperand(1));
+      assert(ArgStructAlloca &&
+             "Unable to find the alloca instruction corresponding to arguments "
+             "for extracted function");
+      StructType *ArgStructType =
+          dyn_cast<StructType>(ArgStructAlloca->getAllocatedType());
+      assert(ArgStructType && "Unable to find struct type corresponding to "
+                              "arguments for extracted function");
+      SharedsSize =
+          Builder.getInt64(M.getDataLayout().getTypeStoreSize(ArgStructType));
+    }
+
+    // Emit the @__kmpc_omp_task_alloc runtime call
+    // The runtime call returns a pointer to an area where the task captured
+    // variables must be copied before the task is run (TaskData)
+    CallInst *TaskData = Builder.CreateCall(
+        TaskAllocFn, {/*loc_ref=*/Ident, /*gtid=*/ThreadID, /*flags=*/Flags,
+                      /*sizeof_task=*/TaskSize, /*sizeof_shared=*/SharedsSize,
+                      /*task_func=*/&OutlinedFn});
+
+    // Get the pointer to loop lb, ub, step from task ptr
+    // and set up the lowerbound,upperbound and step values
+    llvm::Value *lb =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 5);
+    //  Value *LbVal_ext = Builder.CreateSExt(LBVal, Builder.getInt64Ty());
+    Builder.CreateStore(LBVal, lb);
+
+    llvm::Value *ub =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 6);
+    Builder.CreateStore(UBVal, ub);
+
+    llvm::Value *step =
+        Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 7);
+    Value *Step_ext = Builder.CreateSExt(StepVal, Builder.getInt64Ty());
+    Builder.CreateStore(Step_ext, step);
+    llvm::Value *loadstep = Builder.CreateLoad(Builder.getInt64Ty(), step);
+
+    if (HasShareds) {
+      Value *Shareds = StaleCI->getArgOperand(1);
+      Align Alignment = TaskData->getPointerAlignment(M.getDataLayout());
+      Value *TaskShareds = Builder.CreateLoad(VoidPtr, TaskData);
+      Builder.CreateMemCpy(TaskShareds, Alignment, Shareds, Alignment,
+                           SharedsSize);
+    }
+
+    // set up the arguments for emitting kmpc_taskloop runtime call
+    // setting default values for ifval, nogroup, sched, grainsize, task_dup
+    Value *IfVal = Builder.getInt32(1);
+    Value *NoGroup = Builder.getInt32(1);
+    Value *Sched = Builder.getInt32(0);
+    Value *GrainSize = Builder.getInt64(0);
+    Value *TaskDup = Constant::getNullValue(Builder.getPtrTy());
+
+    Value *Args[] = {Ident,    ThreadID, TaskData, IfVal,     lb,     ub,
+                     loadstep, NoGroup,  Sched,    GrainSize, TaskDup};
+
+    // taskloop runtime call
+    Function *TaskloopFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_taskloop);
+    Builder.CreateCall(TaskloopFn, Args);
+
+    // Emit the @__kmpc_end_taskgroup runtime call to end the taskgroup
+    Function *EndTaskgroupFn =
+        getOrCreateRuntimeFunctionPtr(OMPRTL___kmpc_end_taskgroup);
+    Builder.CreateCall(EndTaskgroupFn, {Ident, ThreadID});
+
+    StaleCI->eraseFromParent();
+
+    Builder.SetInsertPoint(TaskloopAllocaBB, TaskloopAllocaBB->begin());
+
+    if (HasShareds) {
+      LoadInst *Shareds = Builder.CreateLoad(VoidPtr, OutlinedFn.getArg(1));
+      OutlinedFn.getArg(1)->replaceUsesWithIf(
+          Shareds, [Shareds](Use &U) { return U.getUser() != Shareds; });
+    }
+
+    Value *IV = CLI->getIndVar();
+    Type *IVTy = IV->getType();
+    Constant *One = ConstantInt::get(IVTy, 1);
+
+    Value *task_lb = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
+                                             OutlinedFn.getArg(1), 5, "gep_lb");
+    Value *LowerBound = Builder.CreateLoad(IVTy, task_lb, "lb");
+
+    Value *task_ub = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
+                                             OutlinedFn.getArg(1), 6, "gep_ub");
+    Value *UpperBound = Builder.CreateLoad(IVTy, task_ub, "ub");
+
+    Builder.SetInsertPoint(CLI->getPreheader()->getTerminator());
+
+    Value *TripCountMinusOne = Builder.CreateSub(UpperBound, LowerBound);
+    Value *TripCount = Builder.CreateAdd(TripCountMinusOne, One, "trip_cnt");
+    // set the trip count in the CLI
+    CLI->setTripCount(TripCount);
+
+    Builder.SetInsertPoint(CLI->getBody(),
+                           CLI->getBody()->getFirstInsertionPt());
+
+    llvm::BasicBlock *Body = CLI->getBody();
+    for (llvm::Instruction &I : *Body) {
+      if (auto *Add = llvm::dyn_cast<llvm::BinaryOperator>(&I)) {
+        if (Add->getOpcode() == llvm::Instruction::Add) {
+          if (llvm::isa<llvm::BinaryOperator>(Add->getOperand(0))) {
+            // update the starting index of the loop
+            Add->setOperand(1, LowerBound);
+          }
+        }
+      }
+    }
+
+    for (Instruction *I : llvm::reverse(ToBeDeleted)) {
+      I->eraseFromParent();
+    }
+  };
+
+  addOutlineInfo(std::move(OI));
+  Builder.SetInsertPoint(TaskloopExitBB, TaskloopExitBB->begin());
+  return Builder.saveIP();
+}
+
 OpenMPIRBuilder::InsertPointOrErrorTy OpenMPIRBuilder::createTask(
     const LocationDescription &Loc, InsertPointTy AllocaIP,
     BodyGenCallbackTy BodyGenCB, bool Tied, Value *Final, Value *IfCondition,
diff --git a/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp b/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
index 8edec990eaaba..d69fcd3db0413 100644
--- a/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
+++ b/mlir/lib/Target/LLVMIR/Dialect/OpenMP/OpenMPToLLVMIRTranslation.cpp
@@ -323,6 +323,18 @@ static LogicalResult checkImplementationStatus(Operation &op) {
     if (op.getDistScheduleChunkSize())
       result = todo("dist_schedule with chunk_size");
   };
+  auto checkFinal = [&todo](auto op, LogicalResult &result) {
+    if (op.getFinal())
+      result = todo("final");
+  };
+  auto checkGrainsize = [&todo](auto op, LogicalResult &result) {
+    if (op.getGrainsize())
+      result = todo("grainsize");
+  };
+  auto checkIf = [](auto op, LogicalResult &) {
+    if (op.getIfExpr())
+      op.emitWarning("if");
+  };
   auto checkHint = [](auto op, LogicalResult &) {
     if (op.getHint())
       op.emitWarning("hint clause discarded");
@@ -340,10 +352,22 @@ static LogicalResult checkImplementationStatus(Operation &op) {
     if (!op.getLinearVars().empty() || !op.getLinearStepVars().empty())
       result = todo("linear");
   };
+  auto checkMergeable = [&todo](auto op, LogicalResult &result) {
+    if (op.getMergeable())
+      result = todo("mergeable");
+  };
+  auto checkNogroup = [&todo](auto op, LogicalResult &result) {
+    if (op.getNogroup())
+      result = todo("nogroup");
+  };
   auto checkNowait = [&todo](auto op, LogicalResult &result) {
     if (op.getNowait())
       result = todo("nowait");
   };
+  auto checkNumTasks = [&todo](auto op, LogicalResult &result) {
+    if (op.getNumTasks())
+      result = todo("num_tasks");
+  };
   auto checkOrder = [&todo](auto op, LogicalResult &result) {
     if (op.getOrder() || op.getOrderMod())
       result = todo("order");
@@ -417,7 +441,15 @@ static LogicalResult checkImplementationStatus(Operation &op) {
         checkNowait(op, result);
       })
       .Case([&](omp::TaskloopOp op) {
-        // TODO: Add other clauses check
+        checkAllocate(op, result);
+        checkFinal(op, result);
+        checkGrainsize(op, result);
+        checkIf(op, result);
+        checkInReduction(op, result);
+        checkMergeable(op, result);
+        checkNogroup(op, result);
+        checkNumTasks(op, result);
+        checkReduction(op, result);
         checkUntied(op, result);
         checkPriority(op, result);
       })
@@ -2097,6 +2129,8 @@ class TaskContextStructManager {
   /// private decls.
   void createGEPsToPrivateVars();
 
+  llvm::Value *isAllocated();
+
   /// De-allocate the task context structure.
   void freeStructPtr();
 
@@ -2177,13 +2211,26 @@ void TaskContextStructManager::createGEPsToPrivateVars() {
   }
 }
 
+llvm::Value *TaskContextStructManager::isAllocated() {
+  if (!structPtr)
+    return nullptr;
+
+  return builder.CreateIsNotNull(structPtr);
+}
+
 void TaskContextStructManager::freeStructPtr() {
   if (!structPtr)
     return;
 
   llvm::IRBuilderBase::InsertPointGuard guard{builder};
-  // Ensure we don't put the call to free() after the terminator
-  builder.SetInsertPoint(builder.GetInsertBlock()->getTerminator());
+  llvm::BasicBlock *currentBlock = builder.GetInsertBlock();
+  if (currentBlock->getTerminator()) {
+    // Ensure we don't put the call to free() after the terminator
+    builder.SetInsertPoint(currentBlock->getTerminator());
+  } else {
+    // Insert the call to free() at the end of the current block
+    builder.SetInsertPoint(currentBlock);
+  }
   builder.CreateFree(structPtr);
 }
 
@@ -2419,6 +2466,207 @@ convertOmpTaskOp(omp::TaskOp taskOp, llvm::IRBuilderBase &builder,
   return success();
 }
 
+// Converts an OpenMP taskloop construct into LLVM IR using OpenMPIRBuilder.
+static LogicalResult
+convertOmpTaskloopOp(Operation &opInst, llvm::IRBuilderBase &builder,
+                     LLVM::ModuleTranslation &moduleTranslation) {
+  using InsertPointTy = llvm::OpenMPIRBuilder::InsertPointTy;
+  auto taskloopOp = cast<omp::TaskloopOp>(opInst);
+  if (failed(checkImplementationStatus(opInst)))
+    return failure();
+
+  // It stores the pointer of allocated firstprivate copies,
+  // which can be used later for freeing the allocated space.
+  SmallVector<llvm::Value *> llvmFirstPrivateVars;
+  PrivateVarsInfo privateVarsInfo(taskloopOp);
+  TaskContextStructManager taskStructMgr{builder, moduleTranslation,
+                                         privateVarsInfo.privatizers};
+
+  llvm::OpenMPIRBuilder::InsertPointTy allocaIP =
+      findAllocaInsertPoint(builder, moduleTranslation);
+
+  assert(builder.GetInsertPoint() == builder.GetInsertBlock()->end());
+  llvm::BasicBlock *taskloopStartBlock = llvm::BasicBlock::Create(
+      builder.getContext(), "omp.taskloop.start",
+      /*Parent=*/builder.GetInsertBlock()->getParent());
+  llvm::Instruction *branchToTaskloopStartBlock =
+      builder.CreateBr(taskloopStartBlock);
+  builder.SetInsertPoint(branchToTaskloopStartBlock);
+
+  llvm::BasicBlock *copyBlock =
+      splitBB(builder, /*CreateBranch=*/true, "omp.private.copy");
+  llvm::BasicBlock *initBlock =
+      splitBB(builder, /*CreateBranch=*/true, "omp.private.init");
+
+  LLVM::ModuleTranslation::SaveStack<OpenMPAllocaStackFrame> frame(
+      moduleTranslation, allocaIP);
+
+  // Allocate and initialize private variables
+  builder.SetInsertPoint(initBlock->getTerminator());
+
+  taskStructMgr.generateTaskContextStruct();
+  taskStructMgr.createGEPsToPrivateVars();
+
+  llvmFirstPrivateVars.resize(privateVarsInfo.blockArgs.size());
+  int index = 0;
+
+  for (auto [privDecl, mlirPrivVar, blockArg, llvmPrivateVarAlloc] :
+       llvm::zip_equal(privateVarsInfo.privatizers, privateVarsInfo.mlirVars,
+                       privateVarsInfo.blockArgs,
+                       taskStructMgr.getLLVMPrivateVarGEPs())) {
+    // To be handled inside the taskloop.
+    if (!privDecl.readsFromMold())
+      continue;
+    assert(llvmPrivateVarAlloc &&
+           "reads from mold so shouldn't have been skipped");
+
+    llvm::Expected<llvm::Value *> privateVarOrErr =
+        initPrivateVar(builder, moduleTranslation, privDecl, mlirPrivVar,
+                       blockArg, llvmPrivateVarAlloc, initBlock);
+    if (!privateVarOrErr)
+      return handleError(privateVarOrErr, *taskloopOp.getOperation());
+
+    llvmFirstPrivateVars[index++] = privateVarOrErr.get();
+
+    llvm::IRBuilderBase::InsertPointGuard guard(builder);
+    builder.SetInsertPoint(builder.GetInsertBlock()->getTerminator());
+
+    if ((privateVarOrErr.get() != llvmPrivateVarAlloc) &&
+        !mlir::isa<LLVM::LLVMPointerType>(blockArg.getType())) {
+      builder.CreateStore(privateVarOrErr.get(), llvmPrivateVarAlloc);
+      // Load it so we have the value pointed to by the GEP
+      llvmPrivateVarAlloc = builder.CreateLoad(privateVarOrErr.get()->getType(),
+                                               llvmPrivateVarAlloc);
+    }
+    assert(llvmPrivateVarAlloc->getType() ==
+           moduleTranslation.convertType(blockArg.getType()));
+  }
+
+  // firstprivate copy region
+  setInsertPointForPossiblyEmptyBlock(builder, copyBlock);
+  if (failed(copyFirstPrivateVars(
+          taskloopOp, builder, moduleTranslation, privateVarsInfo.mlirVars,
+          taskStructMgr.getLLVMPrivateVarGEPs(), privateVarsInfo.privatizers,
+          taskloopOp.getPrivateNeedsBarrier())))
+    return llvm::failure();
+
+  // Set up inserttion point for call to createTaskloop()
+  builder.SetInsertPoint(taskloopStartBlock);
+
+  auto bodyCB = [&](InsertPointTy allocaIP,
+                    InsertPointTy codegenIP) -> llvm::Error {
+    // Save the alloca insertion point on ModuleTranslation stack for use in
+    // nested regions.
+    LLVM::ModuleTranslation::SaveStack<OpenMPAllocaStackFrame> frame(
+        moduleTranslation, allocaIP);
+
+    // translate the body of the taskloop:
+    builder.restoreIP(codegenIP);
+
+    llvm::BasicBlock *privInitBlock = nullptr;
+    privateVarsInfo.llvmVars.resize(privateVarsInfo.blockArgs.size());
+    for (auto [i, zip] : llvm::enumerate(llvm::zip_equal(
+             privateVarsInfo.blockArgs, privateVarsInfo.privatizers,
+             privateVarsInfo.mlirVars))) {
+      auto [blockArg, privDecl, mlirPrivVar] = zip;
+      // This is handled before the task executes
+      if (privDecl.readsFromMold())
+        continue;
+
+      llvm::IRBuilderBase::InsertPointGuard guard(builder);
+      llvm::Type *llvmAllocType =
+          moduleTranslation.convertType(privDecl.getType());
+      builder.SetInsertPoint(allocaIP.getBlock()->getTerminator());
+      llvm::Value *llvmPrivateVar = builder.CreateAlloca(
+          llvmAllocType, /*ArraySize=*/nullptr, "omp.private.alloc");
+
+      llvm::Expected<llvm::Value *> privateVarOrError =
+          initPrivateVar(builder, moduleTranslation, privDecl, mlirPrivVar,
+                         blockArg, llvmPrivateVar, privInitBlock);
+      if (!privateVarOrError)
+        return privateVarOrError.takeError();
+      moduleTranslation.mapValue(blockArg, privateVarOrError.get());
+      privateVarsInfo.llvmVars[i] = privateVarOrError.get();
+      // Add private var to  llvmFirstPrivateVars
+      llvmFirstPrivateVars[index++] = privateVarOrError.get();
+    }
+
+    taskStructMgr.createGEPsToPrivateVars();
+    for (auto [i, llvmPrivVar] :
+         llvm::enumerate(taskStructMgr.getLLVMPrivateVarGEPs())) {
+      if (!llvmPrivVar) {
...
[truncated]

Copy link
Contributor

@tblah tblah left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for posting this Kaviya.

I have a few questions but for the most part I think this should be good to go (with the extra todo message I mention in one of the comments). I understand that you don't have time to work on this immediately so I would suggest merging this without much modification and then improvements can be made in smaller followup patches.

// and set up the lowerbound,upperbound and step values
llvm::Value *lb =
Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop, TaskData, 5);
// Value *LbVal_ext = Builder.CreateSExt(LBVal, Builder.getInt64Ty());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this sign extension needed? Maybe there should just be an assertion here that the types match?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, sign extension is required. I referred Clang’s implementation, where the lower bound, upper bound and step values are defined as kmp_uint64.

 // Build struct kmp_task_t {
  //         void *              shareds;
  //         kmp_routine_entry_t routine;
  //         kmp_int32           part_id;
  //         kmp_cmplrdata_t data1;
  //         kmp_cmplrdata_t data2;
  // For taskloops additional fields:
  //         kmp_uint64          lb;
  //         kmp_uint64          ub;
  //         kmp_int64           st;
  //         kmp_int32           liter;
  //         void *              reductions;
  //       };

// set up the arguments for emitting kmpc_taskloop runtime call
// setting default values for ifval, nogroup, sched, grainsize, task_dup
Value *IfVal = Builder.getInt32(1);
Value *NoGroup = Builder.getInt32(1);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't this default to 0 (aka no clause specified)?

Edit: oh I see we are building the taskgroup explicitly instead of using the runtime. That's okay but I am curious why you decided to do that?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is no specific reason for building the taskgroup explicitly. As mentioned in the above comment, I followed Clang’s implementation, which invokes an explicit taskgroup and aligned the implementation with clang's behaviour.

Comment on lines +2115 to +2122
if (auto *Add = llvm::dyn_cast<llvm::BinaryOperator>(&I)) {
if (Add->getOpcode() == llvm::Instruction::Add) {
if (llvm::isa<llvm::BinaryOperator>(Add->getOperand(0))) {
// update the starting index of the loop
Add->setOperand(1, LowerBound);
}
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why exactly is this needed? I'm worried that this might match other Add instructions?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I agree with your comments that it might match other add instruction. The reason behind doing this change is that the taskloop construct divides the loop iterations into chunks and each chunk is executed as an explicit task. The loop bounds (lower bound, upper bound and step) for these chunks are computed by the runtime function __kmpc_taskloop(...) so we need to update the loop nest with the bounds returned by the runtime.

The current loop-nest translation sets the global loop bounds. This change ensures that the loop bounds are adjusted according to the values returned by the runtime.

I explored several alternative approaches to update the loop-bounds based on runtime, but none of them worked. Also the loop-nest translation is done at this stage where it returns the runtime loop bound values.

Could you share your thoughts, if you have better suggestion for handling this scenario?

};
auto checkIf = [](auto op, LogicalResult &) {
if (op.getIfExpr())
op.emitWarning("if");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this a warning instead of using todo()?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apologies, It should be a TODO check. Let me update it in follow up patch.

Value *NoGroup = Builder.getInt32(1);
Value *Sched = Builder.getInt32(0);
Value *GrainSize = Builder.getInt64(0);
Value *TaskDup = Constant::getNullValue(Builder.getPtrTy());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taskloop works internally by duplicating the current task recursively until there are the right number of tasks to execute the loop with the requested degree of parallelism. I've only skimmed the code but I think it makes a direct copy of the task data structure allocated by the openmp runtime.

I wonder if we will need to use the task dup function for duplicating the task context structure allocated in OpenMPToLLVMIRConversion. Otherwise all of the tasks will share the same pointer to the same task context structure allocated in OpenMPToLLVMIRConversion.

Fixing this will take some work so I suggest we leave it as it is and land this patch emitting TODO when the task context structure isn't empty. Then someone can fix this in a follow up patch.

Does that sound okay to you?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I will add a TODO for it.

// CHECK: br label %taskloop.exit

// CHECK: taskloop.exit:
// CHECK: tail call void @free(ptr %[[OMP_TASK_CONTEXT_PTR]])
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A note to whoever implements nogroup:

The location of this free is valid because end_taskgroup waits until all generated tasks are complete before returning.

If end_taskgroup is not called, some other mechanism will have to be used to ensure that this free is not called until every thread has completed execution.

Copy link
Contributor

@Stylie777 Stylie777 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the patch Kaviya!

I have a few comments but they do not block merging of this patch. Only thing I would like to see is TODO tests for all the TODO messages that have been added. If we need to add those in follow on patch, then thats ok too.


// dummy check to ensure that the task context structure is accessed inside
// the outlined fn.
llvm::Value *cond = taskStructMgr.isAllocated();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unused variable, we can remove this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is dummy check added with purpose.

For taskloop translation, I need to emit a call to __kmpc_taskloop(), which is taking a pointer to __kmpc_omp_task_alloc(). The outlined function _QQmain..omp_par() performs all the computation associated with the task.

Inside _QQmain..omp_par(), %omp.task.context_ptr is checked before extracting the private variables, and this pointer is freed at the end of the function. This logic resulted in a double-free issue with taskloop. To resolve the issue, I moved the deallocation so that %omp.task.context_ptr is freed only after the end of the taskgroup.

However, after moving this logic outside the outlined function, the outlined function _QQmain..omp_par(), no longer receives the second parameter (the pointer to the task structure). So I added a dummy check to check the allocation status of %omp.task.context_ptr, which passes the second parameter in all the cases.

return loopInfo;
};

llvm::OpenMPIRBuilder &ompBuilder = *moduleTranslation.getOpenMPBuilder();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unused variable, we can remove this.

OpenMPIRBuilder::InsertPointOrErrorTy OpenMPIRBuilder::createTaskloop(
const LocationDescription &Loc, InsertPointTy AllocaIP,
BodyGenCallbackTy BodyGenCB,
llvm::function_ref<llvm::Expected<llvm::CanonicalLoopInfo *>()> loopInfo,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The name of loopInfo here does not seem to match that of the header file, this should really be LoopInfo. I also think we could just pass this as a pointer and an assert to check its present, if you disagree let me know.

Suggested change
llvm::function_ref<llvm::Expected<llvm::CanonicalLoopInfo *>()> loopInfo,
llvm::CanonicalLoopInfo * loopInfo,
...
assert(loopInfo && "Expected CLI info");

Type *IVTy = IV->getType();
Constant *One = ConstantInt::get(IVTy, 1);

Value *task_lb = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Naming convention is not consistent with the rest of the file.

Suggested change
Value *task_lb = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
Value *TaskLb = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,

OutlinedFn.getArg(1), 5, "gep_lb");
Value *LowerBound = Builder.CreateLoad(IVTy, task_lb, "lb");

Value *task_ub = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Naming convention is not consistent with rest of file.

Suggested change
Value *task_ub = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,
Value *TaskUb = Builder.CreateStructGEP(OpenMPIRBuilder::Taskloop,


auto loopOp = cast<omp::LoopNestOp>(taskloopOp.getWrappedLoop());

auto loopInfo = [&]() -> llvm::Expected<llvm::CanonicalLoopInfo *> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we could just do the following:

Suggested change
auto loopInfo = [&]() -> llvm::Expected<llvm::CanonicalLoopInfo *> {
llvm::CanonicalLoopInfo *loopInfo{findCurrentLoopInfo(moduleTranslation)};

I am not sure we need a lambda function for this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, it is required because the CLI is available after calling BodyGenCB

 if (Error Err = BodyGenCB(TaskloopAllocaIP, TaskloopBodyIP))
    return Err;

};
auto checkMergeable = [&todo](auto op, LogicalResult &result) {
if (op.getMergeable())
result = todo("mergeable");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to add the TODO tests for all these clauses, as currently only Untied is covered by tests

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, Let me add TODO testcases for all the clauses in a follow-up patch.

@kaviya2510 kaviya2510 marked this pull request as ready for review November 11, 2025 16:57
@kaviya2510
Copy link
Contributor Author

Hi @tblah and @Stylie777, Sorry for the delayed response.
Thankyou for reviewing the patch and sharing your comment. I will start addressing your review comments and update the patch.

@llvmbot llvmbot added flang Flang issues not falling into any other category flang:fir-hlfir labels Nov 16, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

clang:openmp OpenMP related changes to Clang flang:fir-hlfir flang:openmp flang Flang issues not falling into any other category mlir:llvm mlir:openmp mlir

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants